Skip to content

Conversation

@nastra
Copy link
Contributor

@nastra nastra commented Jan 5, 2024

Creating a View exists early in ResolveSessionCatalog, thus this is performing a pre-substitution from CreateView to CreateIcebergView.

Most of the code lives inside CreateV2ViewExec to be aligned with how Spark implements the same functionality for V1 View creation in the CreateViewCommand

@github-actions github-actions bot added the spark label Jan 5, 2024
@nastra nastra added this to the Iceberg 1.5.0 milestone Jan 5, 2024
@nastra nastra force-pushed the spark-view-create-support branch 5 times, most recently from a01dff7 to f128c2a Compare January 8, 2024 22:04
@nastra nastra force-pushed the spark-view-create-support branch 2 times, most recently from c7410ca to ced7c0c Compare January 16, 2024 11:40
plan foreach {
case CreateIcebergView(ResolvedIdentifier(_: ViewCatalog, ident), _, _, _, _, query, _, _) =>
val identifier = Spark3Util.toV1TableIdentifier(ident)
ViewHelper.verifyTemporaryObjectsNotExists(false, identifier, query, Seq.empty)
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ideally we'd have these checks be part of CreateV2ViewExec but by the time we reach CreateV2ViewExec the underlying `query' doesn't contain the info needed to detect these cases here.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the identifier is a ResolvedIdentifier, don't we already know that a temporary view does not exist? That's checked when resolving views. Does the UnresolvedIdentifier has allowTemp set to false?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

what the check does is to make sure that a view doesn't reference a temporary/global view. We particularly test for these 2 cases in createViewReferencingGlobalTempView() / createViewReferencingTempView()

comment: Option[String],
properties: Map[String, String],
originalText: Option[String],
query: LogicalPlan,
Copy link
Contributor

@rdblue rdblue Jan 16, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we want the parsed logical plan here? Is the purpose to run the analyzer rules to make sure that it is valid?

My guess is that this came from the Spark implementation, which probably uses the logical plan here for when there is no SQL text (which would be why originalText is optional). Unless I'm missing something, I think what we should do is not have the parsed query. Instead, I would have a validation that parses and analyzes the query and throws an exception if analysis fails (because of bad references, missing tables, or reliance on temporary views).

}

val queryColumnNames = analyzedPlan.schema.fieldNames
SchemaUtils.checkColumnNameDuplication(queryColumnNames, SQLConf.get.resolver)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this matter if the column aliases were specified?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we want to perform this check, because Spark does the same and fails if you provide duplicate columns in the underlying query:

spark-sql (default)> create temporary view tempv2 as select id, id from iceberg1.foo WHERE id < 12;
[COLUMN_ALREADY_EXISTS] The column `id` already exists. Consider to choose another name or rename the existing column.

I've left that check and also added a test for this

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If there are aliases, we could succeed though.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

After looking into this a bit more, query does correspond to originalText. That's good because we don't have to call the parser separately and this ensures that the query can be analyzed correctly. We should keep this design for the logical plan.

I think there are a few things that we can improve, though. Since the query plan is linked in like this, it will be automatically analyzed so this should already catch things like trying to project a missing column (by the way, we should have a test for that). It will also catch missing relations (tables and other views) but we probably need to handle resolution a bit differently --- we want view resolution to happen just like it would when loading a view. I think that means that we should call our own code from createViewRelation to rewrite identifiers. (Ideally, we could alias here as well, but it may need some special handling if GetColumnByOrdinal doesn't work with an undetermined type.)

Rewriting identifiers needs to happen immediately so that Spark doesn't substitute any temporary views. I think this needs to be done before ResolveViews so we should do it in RewriteViewCommands.

To summarize:

  • RewriteViewCommands should call our code to rewrite identifiers in the parsed query (and alias columns, if possible)
  • If RewriteViewCommands can't apply the aliases, then we will need a rule that applies them once the query is resolved
  • RewriteViewCommands should also check for temporary view references and fail if any are found (and maybe temporary functions as well?)
  • Checking for the right number of columns should be done in the checker (validation phase) if it isn't done already because we use GetColumnByOrdinal for aliases
  • We should drop the query when possible because it shouldn't be needed after the analysis phase. Instead we should get the schema of the query (after aliasing) and pass that along.

@nastra nastra force-pushed the spark-view-create-support branch from ced7c0c to 99a3439 Compare January 17, 2024 14:33

case c@CreateIcebergView(ResolvedIdentifier(_, ident), _, query, columnAliases, columnComments, _, _, _, _, _, _)
if query.resolved && !c.rewritten =>
val rewritten = rewriteIdentifiers(query, ident.asMultipartIdentifier)
Copy link
Contributor

@rdblue rdblue Jan 25, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@nastra, this can't be done here. It needs to be done before any of Spark's rules run. Otherwise, a temporary view may already have been substituted into the plan and could allow the checks to pass even though the view is invalid.

This rule is only responsible for applying the column aliases and comments. Rewriting the identifiers should be done in RewriteViewCommands.


case CreateView(ResolvedView(resolved), userSpecifiedColumns, comment, properties,
Some(queryText), query, allowExisting, replace) =>
verifyTemporaryObjectsDontExist(resolved.identifier, query)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the query needs to have CTE substitution run before running the temporary object check. Otherwise, there could be conflicts between CTE substitution and temporary names. If there is a conflict, the CTE should be used.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

you're absolutely right. I've done that and also added a test

queryText = queryText,
columnAliases = columnAliases,
columnComments = columnComments,
queryColumnNames = queryColumnNames,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think I'd also be fine with not tracking this and passing an empty array through. That's not correct for Spark, but it would work for us since Iceberg doesn't store these.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

technically we don't need to track it in this PR but we'll eventually need it when we show the properties of views and such. I'd probably keep it here but let me know if you'd like me to remove it here and introduce it in an upcoming PR

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It seems reasonable to include it. Let's just leave it as-is since you even have a test for it.

@Test
public void createViewWithCTE() throws NoSuchTableException {
insertRows(10);
String viewName = "simpleViewWithCTE";
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think we want a test for a CTE that conflicts with a temporary table name.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

not sure if you meant a temp view name here (instead of temp table name) but I added a test where a CTE and a temp view both have the same name and when data is being read, the CTE takes precedences

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

That's correct. I did mean temporary view name. Thanks!

case CreateIcebergView(ResolvedIdentifier(_: ViewCatalog, ident), _, query, columnAliases, _,
_, _, _, _, _, _) =>
verifyColumnCount(ident, columnAliases, query)
SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Looks good.

@rdblue rdblue merged commit 3f6e377 into apache:main Jan 26, 2024
@rdblue
Copy link
Contributor

rdblue commented Jan 26, 2024

Merged! Thanks for all the work on this, @nastra! It's great to have this done.

adnanhemani pushed a commit to adnanhemani/iceberg that referenced this pull request Jan 30, 2024
devangjhabakh pushed a commit to cdouglas/iceberg that referenced this pull request Apr 22, 2024
@nastra nastra deleted the spark-view-create-support branch June 3, 2024 12:54
@nastra nastra self-assigned this Nov 25, 2024
@monologuist
Copy link

@nastra Hello, could you please tell me how to correctly create an Iceberg view in Spark?
I followed the instructions on the official website (https://iceberg.apache.org/docs/1.6.1/spark-ddl/#iceberg-views-in-spark) and created a view based on Spark 3.4 and Iceberg 1.6.1. The statement I used is "create view as {query}", and the Iceberg table involved in the "{query}" actually exists. However, there is no metadata information of this view in my HDFS path. It seems that the view I created is a Spark view (I think the correct Iceberg view should be like the example shown here: https://iceberg.apache.org/view-spec/#appendix-a-an-example). I want to create an Iceberg view correctly, but I haven't found more instructions on the official website.
image
image
image

@nastra
Copy link
Contributor Author

nastra commented Apr 10, 2025

@monologuist this looks all correct to me and you should have a JSON file on HDFS that is gzipped and looks similarly to the example from https://iceberg.apache.org/view-spec/#appendix-a-an-example.

@monologuist
Copy link

@monologuist this looks all correct to me and you should have a JSON file on HDFS that is gzipped and looks similarly to the example from https://iceberg.apache.org/view-spec/#appendix-a-an-example.

@nastra Thank you very much for your reply! As you can see, I created a view named my_iceberg_view_3 under the database tpcds_1000_parquet_iceberg_from_text. However, the HDFS path hdfs://xxx/warehouse/tablespace/managed/hive/tpcds_1000_parquet_iceberg_from_text.db/my_iceberg_view_3 does not exist. So this is where I'm confused.

@monologuist
Copy link

@nastra It should be noted that I'm using a Hive catalog. My Spark command is spark-sql --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkSessionCatalog --conf spark.sql.catalog.spark_catalog.type=hive --conf spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions --properties-file ${SPARK_HOME}/conf/spark-defaults.conf --conf spark.sql.catalog.spark_catalog.uri=thrift://xxx:9083. Is the reason I can't create a correct Iceberg view that Iceberg version 1.6.1 doesn't support the view feature for the Hive catalog yet? (#9852)

@nastra
Copy link
Contributor Author

nastra commented Apr 11, 2025

Ah yes, view support for Hive was added in Iceberg 1.7.0

@monologuist
Copy link

monologuist commented Apr 14, 2025

@nastra Thank you for your response! However, when I switched to Iceberg version 1.7.2, the issue remained. Did I make a mistake in my operation?

@nastra
Copy link
Contributor Author

nastra commented Apr 14, 2025

@monologuist can you please open a separate GH issue with detailed repro steps for 1.7.x?

@monologuist
Copy link

@nastra I encounter the error Creating a view is not supported by catalog: spark_catalog when using SparkSessionCatalog.After I switched to Iceberg version 1.7.2, I was able to perform operations related to Iceberg materialized views using the SparkCatalog.
It seems that my problem has been solved for now.

I also want to know that theoretically, Iceberg materialized views support multiple engines([https://iceberg.apache.org/view-spec/#background-and-motivation]), but currently, they are only adapted to Spark, right? I noticed that the official website doesn't mention the support for Iceberg views in Flink, Doris, etc. Is there any plan for when this support will be added?

@nastra
Copy link
Contributor Author

nastra commented Apr 14, 2025

FYI those are not materialized views but normal views. Currently Iceberg view support has been implemented in Spark/Trino. I don't know what's being planned for Flink/Doris or any other engines. Engines like Apache Doris are maintained outside of the Iceberg community, so you will have to check with them what their plan is on supporting Iceberg views.

@monologuist
Copy link

Thank you for your patient explanation. My goal is to make Iceberg views available across multiple engines. I'll check with other communities about their plans in this regard.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

Status: Done

Development

Successfully merging this pull request may close these issues.

3 participants